-
Notifications
You must be signed in to change notification settings - Fork 13.7k
[FLINK-38461] Introduce SinkUpsertMaterializerV2 #27070
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Conversation
This allows to use RowTypeUtils both in planner and in runtime
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @rkhachatryan for the pr!
Overall LGTM, I left some comments, please take a look when you are available, thanks
TABLE_EXEC_SINK_UPSERT_MATERIALIZE_STRATEGY = | ||
key("table.exec.sink.upsert-materialize.strategy") | ||
.enumType(SinkUpsertMaterializeStrategy.class) | ||
.noDefaultValue() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The default is LEGACY
, why noDefaultValue()
here?
"When using strategy=ADAPTIVE, defines the number of entries per key when the implementation is changed from MAP to VALUE. If not specified, Flink uses state-backend specific defaults.") | ||
.linebreak() | ||
.text("The option takes effect during job (re)starting") | ||
.linebreak() | ||
.build()); | ||
|
||
@Documentation.TableOption(execMode = Documentation.ExecMode.STREAMING) | ||
public static final ConfigOption<Long> | ||
TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_HIGH = | ||
key("table.exec.sink.upsert-materialize.adaptive.threshold.high") | ||
.longType() | ||
.noDefaultValue() | ||
.withDescription( | ||
Description.builder() | ||
.text( | ||
"When using strategy=ADAPTIVE, defines the number of entries per key when the implementation is changed from VALUE to MAP. If not specified, Flink uses state-backend specific defaults.") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would you mind mentioning the specific default value for each state backend?
.text( | ||
"When using strategy=ADAPTIVE, defines the number of entries per key when the implementation is changed from VALUE to MAP. If not specified, Flink uses state-backend specific defaults.") | ||
.linebreak() | ||
.text("The option takes effect during job (re)starting") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I do not understand why mentioning "The option takes effect during job (re)starting"
. As I understand, almost all of config options take effect after job (re)starting.
Do you wanna point out that these 2 config options cannot be changed via dynamic conf?
+ Arrays.toString(indexMapping) | ||
+ ", mutableRow=" | ||
+ row | ||
+ '}'; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: how about adding isNullAtNonProjected
into toString()
as well?
import org.apache.flink.table.planner.plan.utils.{AggregateUtil, OverAggregateUtil, SortUtil} | ||
import org.apache.flink.table.planner.plan.utils.PythonUtil.isPythonAggregate | ||
import org.apache.flink.table.planner.utils.ShortcutUtils | ||
import org.apache.flink.table.typeutils.RowTypeUtils |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This file does not change any code, why import is needed? by mistake or do I miss something?
upsertMaterialize, | ||
// persist upsertMaterialize strategy separately in the compiled plan to make it immutable; | ||
// later on, it can't be obtained from the node config because it is merged with the new environment | ||
config.getOptional(TABLE_EXEC_SINK_UPSERT_MATERIALIZE_STRATEGY).orElse(null), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why it can be null?
config.getOptional( | ||
TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_HIGH) | ||
.orElse(null), | ||
config.getOptional( | ||
TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_LOW) | ||
.orElse(null), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
config.getOptional( | |
TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_HIGH) | |
.orElse(null), | |
config.getOptional( | |
TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_LOW) | |
.orElse(null), | |
config.get(TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_HIGH), | |
config.get(TABLE_EXEC_SINK_UPSERT_MATERIALIZE_ADAPTIVE_THRESHOLD_LOW), |
It they are not set, and no default value, get will return null directly.
: HashCodeGenerator.generateRowHash( | ||
new CodeGeneratorContext(config, classLoader), | ||
RowTypeUtils.projectRowType(physicalRowType, inputUpsertKey), | ||
"", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about setting a name here?
TTL_CONFIG, RowType.of(types), EQUALISER, UPSERT_KEY_EQUALISER, upsertKey); | ||
case MAP: | ||
return createV2( | ||
types, | ||
upsertKey, | ||
SequencedMultiSetStateConfig.forMap( | ||
PROCESSING_TIME, StateTtlConfig.DISABLED)); | ||
case VALUE: | ||
return createV2( | ||
types, | ||
upsertKey, | ||
SequencedMultiSetStateConfig.forValue( | ||
PROCESSING_TIME, StateTtlConfig.DISABLED)); | ||
case ADAPTIVE: | ||
return createV2( | ||
types, | ||
upsertKey, | ||
SequencedMultiSetStateConfig.adaptive( | ||
PROCESSING_TIME, 10L, 5L, StateTtlConfig.DISABLED)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
how about extracting a method to create StateTtlConfig
via isTtlSupported()
method?
It is helpful for keep the code consistency, only change isTtlSupported()
in the future while supporting TTL.
? ProjectedRowData.from(inputUpsertKey) | ||
.withNullAtNonProjected(true) | ||
.replaceRow(row) | ||
: row, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have 2 comments about this part since it is a hot code path:
withNullAtNonProjected
will create a newProjectedRowData
object, so it created 2ProjectedRowData
objects for each row.ProjectedRowData
is created on open in SinkUpsertMaterializer[1], and then it is reused by all rows. So how about createProjectedRowData
only once as well here.
[1]
Line 123 in e44d638
upsertKeyProjectedRow2 = ProjectedRowData.from(inputUpsertKey); |
This PR adds a new version of
SinkUpsertMaterializer
based on Sequenced MultiSet state introduced in #27071.The existing operator is not changed.
Test coverage is provided by parameterized the tests extended in #27068.